Skip to content

Conversation

@openinx
Copy link
Member

@openinx openinx commented Sep 4, 2020

This patch will create a separate flink runtime module named flink-runtime, it will shade the common dependency jars and archive all flink connector related classes into a jar. Now I have the basic verification under my localhost as the following:

  1. Downloading the apache flink 1.11 release binary
wget https://www.apache.org/dyn/closer.lua/flink/flink-1.11.1/flink-1.11.1-bin-scala_2.12.tgz
tar xzvf flink-1.11.1-bin-scala_2.12.tgz
cd flink-1.11.1
  1. Start the flink cluster with hadoop environment loaded.
export HADOOP_CLASSPATH=`/Users/openinx/software/hadoop-binary/hadoop-2.9.2/bin/hadoop classpath`
./bin/start-cluster.sh   # start flink cluster
  1. Build the iceberg runtime jar.
<apache-iceberg-project-rootdir>/gradlew build -x test

We will see the runtime jar located like:

 ls -altr /Users/openinx/software/apache-iceberg/flink-runtime/build/libs/iceberg-flink-runtime-850a44c.jar 
-rw-r--r--  1 openinx  staff  34621584 Sep  4 11:24 /Users/openinx/software/apache-iceberg/flink-runtime/build/libs/iceberg-flink-runtime-850a44c.jar
  1. Start the flink sql client
# Switch to flink binary root dir

export HADOOP_CLASSPATH=`/Users/openinx/software/hadoop-binary/hadoop-2.9.2/bin/hadoop classpath`
./bin/sql-client.sh \
    embedded \
    -j /Users/openinx/software/apache-iceberg/flink-runtime/build/libs/iceberg-flink-runtime-850a44c.jar \
    shell
  1. Let's execute few flink sql
Flink SQL> create catalog iceberg_catalog with(
>   'type'='iceberg',
>   'catalog-type'='hadoop',
>   'property-version'='1',
>   'warehouse'='/Users/openinx/software/flink/build-target/hadoop-warehouse'
> );
[INFO] Catalog has been created.

Flink SQL> USE catalog iceberg_catalog;

Flink SQL> CREATE TABLE test (
>     id bigint,
>     data string
> );
[INFO] Table has been created.

Flink SQL> insert into test select 1, 'hello';
[INFO] Submitting SQL update statement to the cluster...
[INFO] Table update statement has been successfully submitted to the cluster:
Job ID: 458cf238116135db262ec7dbec47f32e
  1. Check the iceberg table:
➜  default git:(master) ✗ pwd     
/Users/openinx/software/flink/build-target/hadoop-warehouse/default
➜  default git:(master) ✗ tree -a
.
└── test
    ├── data
    │   ├── .00000-0-bcfe3440-c326-4d17-be1f-ca3056a45376-00001.parquet.crc
    │   └── 00000-0-bcfe3440-c326-4d17-be1f-ca3056a45376-00001.parquet
    └── metadata
        ├── .2220c21a-38ee-4c27-a498-521de74f7eb7-m0.avro.crc
        ├── .snap-8091339106931514070-1-2220c21a-38ee-4c27-a498-521de74f7eb7.avro.crc
        ├── .v1.metadata.json.crc
        ├── .v2.metadata.json.crc
        ├── .version-hint.text.crc
        ├── 2220c21a-38ee-4c27-a498-521de74f7eb7-m0.avro
        ├── snap-8091339106931514070-1-2220c21a-38ee-4c27-a498-521de74f7eb7.avro
        ├── v1.metadata.json
        ├── v2.metadata.json
        └── version-hint.text

3 directories, 12 files

It will need still more work to fill the LICENSE and NOTICE in flink-runtime module, and I will test more cases to confirm whether it works well.

@openinx openinx marked this pull request as draft September 4, 2020 04:14
@openinx
Copy link
Member Author

openinx commented Sep 4, 2020

I used this patch to copy all the dependencies from flink module to a separate folder.

diff --git a/build.gradle b/build.gradle
index 2ee676de..d0d21ec9 100644
--- a/build.gradle
+++ b/build.gradle
@@ -297,6 +297,11 @@ project(':iceberg-flink') {
       exclude group: 'com.zaxxer', module: 'HikariCP'
     }
   }
+
+  task copyDependencies(type: Copy) {
+    from configurations.default
+    into 'dependencies'
+  }
 }
 
 project(':iceberg-flink-runtime') {

Here are the dependencies:

➜  dependencies git:(flink-runtime) ✗ ls -atlr  
total 72120
drwxr-xr-x   6 openinx  staff       192 Sep  4 17:35 ..
-rw-r--r--   1 openinx  staff     70648 Sep  4 17:35 iceberg-data-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff     80259 Sep  4 17:35 iceberg-orc-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff    269140 Sep  4 17:35 iceberg-parquet-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff     31033 Sep  4 17:35 iceberg-hive-metastore-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff    723066 Sep  4 17:35 iceberg-core-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff    341213 Sep  4 17:35 iceberg-api-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff     31339 Sep  4 17:35 iceberg-common-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff    587956 Sep  4 17:35 avro-1.9.2.jar
-rw-r--r--   1 openinx  staff   1781264 Sep  4 17:35 orc-core-1.6.3-nohive.jar
-rw-r--r--   1 openinx  staff     28953 Sep  4 17:35 orc-shims-1.6.3.jar
-rw-r--r--   1 openinx  staff  19512529 Sep  4 17:35 parquet-avro-1.11.0.jar
-rw-r--r--   1 openinx  staff    437637 Sep  4 17:35 parquet-hadoop-1.11.0.jar
-rw-r--r--   1 openinx  staff   1619439 Sep  4 17:35 parquet-column-1.11.0.jar
-rw-r--r--   1 openinx  staff    848718 Sep  4 17:35 parquet-encoding-1.11.0.jar
-rw-r--r--   1 openinx  staff     96197 Sep  4 17:35 parquet-common-1.11.0.jar
-rw-r--r--   1 openinx  staff    700667 Sep  4 17:35 parquet-format-structures-1.11.0.jar
-rw-r--r--   1 openinx  staff     41203 Sep  4 17:35 slf4j-api-1.7.25.jar
-rw-r--r--   1 openinx  staff     15322 Sep  4 17:35 findbugs-annotations-1.3.9-1.jar
-rw-r--r--   1 openinx  staff   1634977 Sep  4 17:35 iceberg-bundled-guava-746c0c8.dirty.jar
-rw-r--r--   1 openinx  staff   1403755 Sep  4 17:35 jackson-databind-2.10.2.jar
-rw-r--r--   1 openinx  staff    349074 Sep  4 17:35 jackson-core-2.10.2.jar
-rw-r--r--   1 openinx  staff    846706 Sep  4 17:35 caffeine-2.7.0.jar
-rw-r--r--   1 openinx  staff    615064 Sep  4 17:35 commons-compress-1.19.jar
-rw-r--r--   1 openinx  staff    193908 Sep  4 17:35 checker-qual-2.6.0.jar
-rw-r--r--   1 openinx  staff     13161 Sep  4 17:35 error_prone_annotations-2.3.3.jar
-rw-r--r--   1 openinx  staff    181098 Sep  4 17:35 aircompressor-0.15.jar
-rw-r--r--   1 openinx  staff    102244 Sep  4 17:35 jaxb-api-2.2.11.jar
-rw-r--r--   1 openinx  staff     19014 Sep  4 17:35 annotations-17.0.0.jar
-rw-r--r--   1 openinx  staff    233745 Sep  4 17:35 threeten-extra-1.5.0.jar
-rw-r--r--   1 openinx  staff     68080 Sep  4 17:35 jackson-annotations-2.10.2.jar
-rw-r--r--   1 openinx  staff   1786250 Sep  4 17:35 parquet-jackson-1.11.0.jar
-rw-r--r--   1 openinx  staff   2021167 Sep  4 17:35 snappy-java-1.1.7.3.jar
-rw-r--r--   1 openinx  staff    111119 Sep  4 17:35 commons-pool-1.6.jar
-rw-r--r--   1 openinx  staff     26586 Sep  4 17:35 javax.annotation-api-1.3.2.jar
drwxr-xr-x  37 openinx  staff      1184 Sep  4 17:35 .
-rw-r--r--   1 openinx  staff     20924 Sep  4 17:35 audience-annotations-0.11.0.jar

@chenjunjiedada
Copy link
Collaborator

Do we expect the Flink cluster has hive execution environment like Spark cluster? In our Flink cluster, we don't have a hive environment so that we require users to upload the application jar along with a hive-exec jar. The spark cluster doesn't need this since it has built-in hive jars. Thoughts?

@openinx
Copy link
Member Author

openinx commented Sep 7, 2020

I'm not quite sure whether we need to pack the hive-exec jar within the flink-runtime jar now. Seems it's a fat jar and include an old version guava dependency ( just notice the comment in build.gradle), I will try the demo under hive environment.

@openinx
Copy link
Member Author

openinx commented Sep 8, 2020

I read the apache flink hive document here, It's recommended to use the bundled flink-sql-connector-hive-x.x.x jar in flink sql client if the hive version is included in bundled list, otherwise we could choose the jars from hive by ourself.

So I think we don't need to include the hive-exec jar within the flink-runtime module again, FYI @chenjunjiedada .

@chenjunjiedada
Copy link
Collaborator

Thanks @openinx! I wasn't aware Flink has this bundle jar.

@rdblue
Copy link
Contributor

rdblue commented Sep 9, 2020

@kbendick, should this be labelled build? The label isn't there, so maybe we're missing build.gradle in the config?

@kbendick
Copy link
Contributor

kbendick commented Sep 13, 2020

@kbendick, should this be labelled build? The label isn't there, so maybe we're missing build.gradle in the config?

Hmm yes it definitely should be. I have **gradle** under BUILD, but I realize now that it should be *gradle*. I will open an issue and then submit a PR. I'm also opening an issue and PR for proactively adding flink-runtime to the flink label @rdblue

@rdblue rdblue merged commit bd37a47 into apache:master Sep 16, 2020
@rdblue
Copy link
Contributor

rdblue commented Sep 16, 2020

Thanks, @openinx! Looks good now.

@rdblue rdblue added this to the Java 0.10.0 Release milestone Nov 16, 2020
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants